518789
@@ -35,6 +35,7 @@
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.locks.ReentrantLock;
 
+import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.common.io.encoded.MemoryBuffer;
 import org.apache.hadoop.hive.conf.HiveConf;
@@ -335,23 +336,28 @@
public long getMaxCacheSize() {
 
   private ByteBuffer preallocate(int arenaSize) {
     if (isMapped) {
+      RandomAccessFile rwf = null;
+      File rf = null;
       Preconditions.checkArgument(isDirect, "All memory mapped allocations have to be direct buffers");
       try {
-        File rf = File.createTempFile("arena-", ".cache", cacheDir.toFile());
-        RandomAccessFile rwf = new RandomAccessFile(rf, "rw");
+        rf = File.createTempFile("arena-", ".cache", cacheDir.toFile());
+        rwf = new RandomAccessFile(rf, "rw");
         rwf.setLength(arenaSize); // truncate (TODO: posix_fallocate?)
         // Use RW, not PRIVATE because the copy-on-write is irrelevant for a deleted file
         // see discussion in YARN-5551 for the memory accounting discussion
         ByteBuffer rwbuf = rwf.getChannel().map(MapMode.READ_WRITE, 0, arenaSize);
-        // A mapping, once established, is not dependent upon the file channel that was used to
-        // create it. delete file and hold onto the map
-        rwf.close();
-        rf.delete();
         return rwbuf;
       } catch (IOException ioe) {
         LlapIoImpl.LOG.warn("Failed trying to allocate memory mapped arena", ioe);
         // fail similarly when memory allocations fail
         throw new OutOfMemoryError("Failed trying to allocate memory mapped arena: " + ioe.getMessage());
+      } finally {
+        // A mapping, once established, is not dependent upon the file channel that was used to
+        // create it. delete file and hold onto the map
+        IOUtils.closeQuietly(rwf);
+        if (rf != null) {
+          rf.delete();
+        }
       }
     }
     return isDirect ? ByteBuffer.allocateDirect(arenaSize) : ByteBuffer.allocate(arenaSize);
